KAFKA-14478: Move LogConfig/CleanerConfig and related to storage module#13049
KAFKA-14478: Move LogConfig/CleanerConfig and related to storage module#13049ijuma merged 7 commits intoapache:trunkfrom
Conversation
9dbdb87 to
9532d07
Compare
There was a problem hiding this comment.
Removed the trim from here since we do it in ensureValid as well.
aaed51e to
4277ed7
Compare
…MetadataLog` (#13051) `LogConfig.validateValues` may fail or incorrectly succeed if the properties don't include defaults. During the conversion of `LogConfig` to Java (#13049), it became clear that the `asInstanceOf[Long]` calls in `LogConfig.validateValues` were converting `null` to `0` when this method was invoked from `KafkaMetadataLog`. This means that it would be possible for it to validate successfully in cases where it should not. Reviewers: José Armando García Sancio <jsancio@apache.org>
6bf3b84 to
f5946bd
Compare
| maxBatchSizeInBytes: Int, | ||
| maxFetchSizeInBytes: Int, | ||
| fileDeleteDelayMs: Int, | ||
| fileDeleteDelayMs: Long, |
There was a problem hiding this comment.
Changed to long to match the type of the config (even if the default value happened to fit an int before).
| .define(LogIndexIntervalBytesProp, INT, LogConfig.DEFAULT_INDEX_INTERVAL_BYTES, atLeast(0), MEDIUM, LogIndexIntervalBytesDoc) | ||
| .define(LogFlushIntervalMessagesProp, LONG, LogConfig.DEFAULT_FLUSH_MESSAGES_INTERVAL, atLeast(1), HIGH, LogFlushIntervalMessagesDoc) | ||
| .define(LogDeleteDelayMsProp, LONG, LogConfig.DEFAULT_FILE_DELETE_DELAY_MS, atLeast(0), HIGH, LogDeleteDelayMsDoc) | ||
| .define(LogFlushSchedulerIntervalMsProp, LONG, LogConfig.DEFAULT_FLUSH_MS, HIGH, LogFlushSchedulerIntervalMsDoc) |
There was a problem hiding this comment.
It's a bit unintuitive that LogFlushSchedulerIntervalMsProp (vs LogFlushIntervalMsProp) defaults to LogConfig.DEFAULT_FLUSH_MS, but this preserves the existing approach. There is logic to set LogFlushIntervalMsProp to LogFlushSchedulerIntervalMsProp if unset.
|
JDK 8 and JDK 11 builds passed, JDK 17 had one unrelated failure. |
| if (segmentJitterMs == 0) | ||
| return 0; | ||
| else | ||
| return Utils.abs(ThreadLocalRandom.current().nextInt()) % Math.min(segmentJitterMs, segmentMs); |
There was a problem hiding this comment.
Slight change here: we now use a ThreadLocalRandom instead of a global Random.
| TopicConfig.LOCAL_LOG_RETENTION_MS_CONFIG, | ||
| TopicConfig.LOCAL_LOG_RETENTION_BYTES_CONFIG, | ||
| LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG, | ||
| FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG |
There was a problem hiding this comment.
Added LEADER_REPLICATION_THROTTLED_REPLICAS_CONFIG and FOLLOWER_REPLICATION_THROTTLED_REPLICAS_CONFIG to this list since they never had server defaults.
| } | ||
|
|
||
| // Visible for testing | ||
| public static class LogConfigDef extends ConfigDef { |
There was a problem hiding this comment.
Removed define overrides since we now rely on the statically defined ServerTopicConfigSynonyms to retrieve the server config default/synonym.
|
@satishd If you have cycles, this PR is ready for review. |
mimaison
left a comment
There was a problem hiding this comment.
Thanks. I left a few small suggestions.
There was a problem hiding this comment.
Maybe put it with the other java.util imports below
| "all replicas for this topic."; | ||
|
|
||
| @SuppressWarnings("deprecation") | ||
| private static final String MESSAGE_FORMAT_VERSION_DOC = TopicConfig.MESSAGE_FORMAT_VERSION_DOC; |
There was a problem hiding this comment.
Do we need this constant? We already suppress deprecation in the static block below.
There was a problem hiding this comment.
We do, there's no way to suppress deprecations within the static block (I made this change to get the code to compile). If you find a way to avoid this constant, please do share.
| private final int maxMessageSize; | ||
| private final Map<?, ?> props; | ||
|
|
||
| public LogConfig(Map<?, ?> props) { |
There was a problem hiding this comment.
Considering the number of callers with new Properties() in tests I wonder if we should have a zero argument constructor too.
There was a problem hiding this comment.
I would prefer to leave it as is. Generally, I don't think it's an improvement overall to add overloads that are only used in tests.
|
@mimaison Thanks for the review. I addressed or replied to each comment. |
| validateValues(valueMaps); | ||
| } | ||
|
|
||
| public static void main(String[] args) { |
There was a problem hiding this comment.
Since we moved the package of this tool, should we change the following line in build.gradle?
mainClass = 'kafka.log.LogConfig'
There was a problem hiding this comment.
I observed this issue when I pulled the trunk changes in other PR and raised a fix for that in #13079
| return Utils.mkEntry(topicConfigName, synonymsWithPrefix); | ||
| } | ||
|
|
||
| private static Entry<String, List<ConfigSynonym>> single(String topicConfigName, String brokerConfigName) { |
There was a problem hiding this comment.
Would differentName be more intuitive than single?
There was a problem hiding this comment.
Good question. The naming scheme was trying to cover 3 different cases, each with prefix/no prefix variants:
- Single mapping between topic and broker config name:
single(this is likeCollections.singletonin a way) - Mapping of topic to multiple broker synonyms:
list - A mapping where the topic and broker config have the same name:
sameName(this could also have been calledidentitynow that I think of it).
Are you suggesting that all single* methods become differentName*? What about list*, does that become differentNameList*?
There was a problem hiding this comment.
Thanks for the explanation. Got it. I am not sure that I could make the naming better. We can just keep it as it is.
…MetadataLog` (apache#13051) `LogConfig.validateValues` may fail or incorrectly succeed if the properties don't include defaults. During the conversion of `LogConfig` to Java (apache#13049), it became clear that the `asInstanceOf[Long]` calls in `LogConfig.validateValues` were converting `null` to `0` when this method was invoked from `KafkaMetadataLog`. This means that it would be possible for it to validate successfully in cases where it should not. Reviewers: José Armando García Sancio <jsancio@apache.org>
…le (apache#13049) Additional notable changes to fix multiple dependency ordering issues: * Moved `ConfigSynonym` to `server-common` * Moved synonyms from `LogConfig` to `ServerTopicConfigSynonyms ` * Removed `LogConfigDef` `define` overrides and rely on `ServerTopicConfigSynonyms` instead. * Moved `LogConfig.extractLogConfigMap` to `KafkaConfig` * Consolidated relevant defaults from `KafkaConfig`/`LogConfig` in the latter * Consolidate relevant config name definitions in `TopicConfig` * Move `ThrottledReplicaListValidator` to `storage` Reviewers: Satish Duggana <satishd@apache.org>, Mickael Maison <mickael.maison@gmail.com>
Additional notable changes to fix multiple dependency ordering issues:
ConfigSynonymtoserver-commonLogConfigtoServerTopicConfigSynonymsLogConfigDefdefineoverrides and rely onServerTopicConfigSynonymsinstead.LogConfig.extractLogConfigMaptoKafkaConfigKafkaConfig/LogConfigin the latterTopicConfigThrottledReplicaListValidatortostorageCommitter Checklist (excluded from commit message)